Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: support COUNT() #11229

Merged
merged 14 commits into from
Jul 17, 2024
Merged

feat: support COUNT() #11229

merged 14 commits into from
Jul 17, 2024

Conversation

tshauck
Copy link
Contributor

@tshauck tshauck commented Jul 2, 2024

Which issue does this PR close?

Closes #11228

Rationale for this change

Adds an analyzer rule that roughly matches the current CountWildcardRule rule. I made them separate, but at first blush they could be combined.

I'm not sure how best to handle expressions in rust-land however, because right now the count function takes an expression as an input.

What changes are included in this PR?

  • adds CountEmptyRule
  • basic slt tests
  • quick cleanup on CountWildcardRule that this is modeled after

Are these changes tested?

Added some slt tests, though need to add more.

Are there any user-facing changes?

@github-actions github-actions bot added optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt) labels Jul 2, 2024

/// Rewrite `Count()` to `Count(Expr:Literal(1))`.
#[derive(Default)]
pub struct CountEmptyRule {}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is heavily inspired/copied from CountWildcardRule. It should be possible to also combine this with that if that's a better path.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As each optimizer pass does have non trivial overhead (like it walks the plan trees) I think it would be better if this was combined into the existing CountWildcardRule if possible

@tshauck tshauck marked this pull request as ready for review July 3, 2024 00:29
@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 3, 2024

I can think of two alternative approaches

  1. We can just add wildcard to count if we found that args is empty
select_exprs: [AggregateFunction(AggregateFunction { func_def: UDF(AggregateUDF { inner: Count { name: "count", signature: Signature { type_signature: VariadicAny, volatility: Immutable } } }), args: [Wildcard { qualifier: None }], distinct: false, filter: None, order_by: None, null_treatment: None })]

select_exprs: [AggregateFunction(AggregateFunction { func_def: UDF(AggregateUDF { inner: Count { name: "count", signature: Signature { type_signature: VariadicAny, volatility: Immutable } } }), args: [], distinct: false, filter: None, order_by: None, null_treatment: None })]

let select_exprs = self.prepare_select_exprs(

  1. We could probably do even more, we convert args to const 1 if there is wildcard or is empty, so we don't even need the optimize rule count_wildcard_rule

I think we could rely on the planner trait we recently added in #11180 , so we can rewrite it with expected args based on the expression we got in sql parser.

@tshauck What do you think?

@tshauck
Copy link
Contributor Author

tshauck commented Jul 3, 2024

@jayzhan211 thanks for the feedback. I like the idea because it would seem to avoid having to update the count function to accept zero args. The current approach is nice at least in that it's very clear what's happening (at least to me) and while a bit of code it's mostly a single chunk.

Can you say more about new planner trait and how that'd be used? Is the idea I'd implement UserDefinedSQLPlanner for a new struct like CountPlanner?

Also, since this is somewhat inspired from duckdb, they do a count_star() function that is distinct from other counts and is an alias for count(). This approach may be nice in that it'd afford the opportunity to have a dataframe expression itself and also wouldn't require updating the args for count.

@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 4, 2024

@jayzhan211 thanks for the feedback. I like the idea because it would seem to avoid having to update the count function to accept zero args. The current approach is nice at least in that it's very clear what's happening (at least to me) and while a bit of code it's mostly a single chunk.

Can you say more about new planner trait and how that'd be used? Is the idea I'd implement UserDefinedSQLPlanner for a new struct like CountPlanner?

Also, since this is somewhat inspired from duckdb, they do a count_star() function that is distinct from other counts and is an alias for count(). This approach may be nice in that it'd afford the opportunity to have a dataframe expression itself and also wouldn't require updating the args for count.

I think we can have a more general Planner, AggregateFunctionPlanner

            if let Some(fm) = self.context_provider.get_aggregate_meta(&name) {
                let order_by = self.order_by_to_sort_expr(
                    &order_by,
                    schema,
                    planner_context,
                    true,
                    None,
                )?;
                let order_by = (!order_by.is_empty()).then_some(order_by);
                let args = self.function_args_to_expr(args, schema, planner_context)?;
                let filter: Option<Box<Expr>> = filter
                    .map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context))
                    .transpose()?
                    .map(Box::new);

                let raw_aggregate_function = RawAggregateFunction {
                    fm,
                    args,
                    distinct,
                    filter,
                    order_by,
                    null_treatment,
                }
                // general planner for rewriting aggregate function
                // we convert  wildcard and empty args to lit(1) for count in our case
                plan_aggregate_function(raw_aggregate_function)

                return Ok(Expr::AggregateFunction(expr::AggregateFunction::new_udf(
                    fm,
                    args,
                    distinct,
                    filter,
                    order_by,
                    null_treatment,
                )));
            }

// User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function
if let Some(fm) = self.context_provider.get_aggregate_meta(&name) {
let order_by = self.order_by_to_sort_expr(
&order_by,
schema,
planner_context,
true,
None,
)?;
let order_by = (!order_by.is_empty()).then_some(order_by);
let args = self.function_args_to_expr(args, schema, planner_context)?;
let filter: Option<Box<Expr>> = filter
.map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context))
.transpose()?
.map(Box::new);
return Ok(Expr::AggregateFunction(expr::AggregateFunction::new_udf(
fm,
args,
distinct,
filter,
order_by,
null_treatment,
)));

This approach may be nice in that it'd afford the opportunity to have a dataframe expression itself and also wouldn't require updating the args for count

I'm not sure which approach are you mentioning and what do you mean about dataframe expression 😕 Are you suggesting we support count_start() dataframe API?

I didn't find count_star() in duckdb, so not sure should we support this
https://duckdb.org/docs/sql/expressions/star.html

we have dataframe API count() and count_distinct() now. If we need count(*), we can use count() with wildcard argument.

@tshauck
Copy link
Contributor Author

tshauck commented Jul 4, 2024

Thanks, your original reply was a bit confusing, but I think that makes sense. Adding something new/generic was more than I had originally had in mind, but I'll take a look at adding something like you mention in AggregateFunctionPlanner.

Here's what I meant w.r.t. duckdb...

D EXPLAIN SELECT COUNT(*)
  ;

┌─────────────────────────────┐
│┌───────────────────────────┐│
││       Physical Plan       ││
│└───────────────────────────┘│
└─────────────────────────────┘
┌───────────────────────────┐
│    UNGROUPED_AGGREGATE    │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│        count_star()       │
└─────────────┬─────────────┘                             
┌─────────────┴─────────────┐
│         DUMMY_SCAN        │
└───────────────────────────┘

You're right on the dataframe you can do count with wildcard, but it doesn't seem as analogous to COUNT().

I'm traveling through the end of the weekend, but'll follow up when I've had a chance to try this out.

@jayzhan211 jayzhan211 mentioned this pull request Jul 5, 2024
@tshauck tshauck force-pushed the support-count-with-no-args branch from cba60ac to 07bee49 Compare July 5, 2024 20:58
@github-actions github-actions bot added sql SQL Planner logical-expr Logical plan and expressions core Core DataFusion crate and removed optimizer Optimizer rules labels Jul 5, 2024
@tshauck
Copy link
Contributor Author

tshauck commented Jul 5, 2024

@jayzhan211 -- would you mind taking a quick look now to see if this approach is inline w/ what you were thinking? It's super rough, but just hoping to get feedback on the general shape.

On this branch I can do:

arrow-datafusion/datafusion-cli support-… ➜ cargo run                                                  
    Finished `dev` profile [unoptimized + debuginfo] target(s) in 0.25s
     Running `target/debug/datafusion-cli`
DataFusion CLI v39.0.0
> SELECT COUNT(*) AS cnt;
+-----+
| cnt |
+-----+
| 1   |
+-----+
1 row(s) fetched. 
Elapsed 0.025 seconds.

> SELECT COUNT() AS cnt;
+-----+
| cnt |
+-----+
| 1   |
+-----+
1 row(s) fetched. 
Elapsed 0.005 seconds.

@jayzhan211
Copy link
Contributor

@jayzhan211 -- would you mind taking a quick look now to see if this approach is inline w/ what you were thinking? It's super rough, but just hoping to get feedback on the general shape.

On this branch I can do:

arrow-datafusion/datafusion-cli support-… ➜ cargo run                                                  
    Finished `dev` profile [unoptimized + debuginfo] target(s) in 0.25s
     Running `target/debug/datafusion-cli`
DataFusion CLI v39.0.0
> SELECT COUNT(*) AS cnt;
+-----+
| cnt |
+-----+
| 1   |
+-----+
1 row(s) fetched. 
Elapsed 0.025 seconds.

> SELECT COUNT() AS cnt;
+-----+
| cnt |
+-----+
| 1   |
+-----+
1 row(s) fetched. 
Elapsed 0.005 seconds.

It is what I'm thinking about!

@tshauck tshauck force-pushed the support-count-with-no-args branch from 6782364 to 4550dbd Compare July 7, 2024 15:49
return Ok(PlannerResult::Planned(Expr::AggregateFunction(
expr::AggregateFunction::new_udf(
aggregate_function.udf,
vec![lit(COUNT_STAR_EXPANSION).alias("*")],
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Quickly tried a few options for aliasing, but nothing quite works right. Need to look a little deeper unless it's obvious to someone else.

@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 10, 2024

Can you say more about how this would work? If this adds complexity, it's not clear to me it's worth it vs the rule. Like why is this better than planning via a logical optimizer rule?

I think the idea is to reduce the complexity, count(), count(1), and count(*) are three equivalent things. Instead of relying on optimizer rules to convert these variations, we could standardize them to a single form, such as count(1), during the query planning stage

Not sure what exactly is count_star you mentioned

IMO, introducing a new count_start function is unnecessary. We already have the capability to handle count(1) in our AggregateStatistics module. Instead, our focus should be on converting count() and count(*) to count(1).

Can you say more about how this would work?

If you are asking dataframe API specifically, it should be something like

pub fn count(expr: Expr) -> Expr {
    let expr = if expr == Expr::Wildcard {
        expr = lit(1)
    } else {
        expr
    }

    Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf(
        count_udaf(),
        vec![expr],
        false,
        None,
        None,
        None,
    ))
}

@tshauck
Copy link
Contributor Author

tshauck commented Jul 10, 2024

I think we both agree on the outcome, I'm talking about the implementation and particular how it works for the alias issues I saw. E.g. your example would break a lot of tests though it'd produce the right answer because it changes how the plan is represented. So the additional complexity I'm talking about is what changes need to support that (if any) and the extent to which that makes it a more complex issue than something like count star which may feel verbose, but seems simple.

This is also an interesting test, because it goes straight to the logical plan builder...

let sql_results = ctx
    .sql("select count(*) from t1")
    .await?
    .select(vec![col("count(*)")])?
    .explain(false, false)?
    .collect()
    .await?;

which maybe I missed it, but wouldn't include potentially being planned?

@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 10, 2024

I think we both agree on the outcome, I'm talking about the implementation and particular how it works for the alias issues I saw. E.g. your example would break a lot of tests though it'd produce the right answer because it changes how the plan is represented. So the additional complexity I'm talking about is what changes need to support that (if any) and the extent to which that makes it a more complex issue than something like count star which may feel verbose, but seems simple.

This is also an interesting test, because it goes straight to the logical plan builder...

let sql_results = ctx
    .sql("select count(*) from t1")
    .await?
    .select(vec![col("count(*)")])?
    .explain(false, false)?
    .collect()
    .await?;

which maybe I missed it, but wouldn't include potentially being planned?

.select(vec![col("count(*)")])?

It seems like a duplicated select. Is this reasonable 🤔

If the alias issue is what I think of, we can add display_name() in AggregateUDFImpl so we can get the name count_star. The idea is similar to display_name in ScalarUDFImpl.

Expr::ScalarFunction(fun) => {
w.write_str(fun.func.display_name(&fun.args)?.as_str())?;
}

break a lot of tests

If only the function displayed name is changed, I think it is acceptable.

@tshauck
Copy link
Contributor Author

tshauck commented Jul 10, 2024

The test is from here:

async fn test_count_wildcard_on_aggregate() -> Result<()> {
let ctx = create_join_context()?;
register_alltypes_tiny_pages_parquet(&ctx).await?;
let sql_results = ctx
.sql("select count(*) from t1")
.await?
.select(vec![col("count(*)")])?
.explain(false, false)?
.collect()
.await?;
// add `.select(vec![count(wildcard())])?` to make sure we can analyze all node instead of just top node.
let df_results = ctx
.table("t1")
.await?
.aggregate(vec![], vec![count(wildcard())])?
.select(vec![count(wildcard())])?
.explain(false, false)?
.collect()
.await?;
//make sure sql plan same with df plan
assert_eq!(
pretty_format_batches(&sql_results)?.to_string(),
pretty_format_batches(&df_results)?.to_string()
);
Ok(())
}
, and while funky, the important part is select(col("count(*)")) which again seems to go directly to the logical planner builder and thus isn't an expression to be planned, or is that right?

@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 10, 2024

the important part is select(col("count(*)")) which again seems to go directly to the logical planner builder and thus isn't an expression to be planned, or is that right?

I agree with it. And, I'm not sure why the test looks like this and whether we should consider it valid or not.

IMO, .select(vec![count(wildcard())])? is the correct way to build the same plan.

I play around it and have a workaround solution for aliasing issue, I guess we can utilize display_name to further simplify the cast and alias

https://github.com/tshauck/arrow-datafusion/pull/1/files#diff-4a599584dfc900ec21169f4f820a1b1db46b004b77533dab83a6178d5d3a467e

If we need to deal with select(col("count(*)")), then we can't just manipulate the expr in ExprPlanner and dataframe API but optimizer. 🤔 And, it also means most of the ExprPlanner we have now are not applied to this kind of plan builder too.

@jayzhan211
Copy link
Contributor

jayzhan211 commented Jul 12, 2024

I found the function name issue is a lot complex than I thought. I think we should have a different name for function for displayed and planning.

In DuckDB, count_star is used across the plan, but

D select count(1) from t;
┌──────────┐
│ count(1) │
│  int64   │
├──────────┤
│        0 │
└──────────┘
D explain select count(1) from t;

┌─────────────────────────────┐
│┌───────────────────────────┐│
││       Physical Plan       ││
│└───────────────────────────┘│
└─────────────────────────────┘
┌───────────────────────────┐
│    UNGROUPED_AGGREGATE    │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│        count_star()       │
└─────────────┬─────────────┘                             
┌─────────────┴─────────────┐
│         SEQ_SCAN          │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│             t             │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│           EC: 0           │
└───────────────────────────┘                             
D
D explain select count(*) from t;

┌─────────────────────────────┐
│┌───────────────────────────┐│
││       Physical Plan       ││
│└───────────────────────────┘│
└─────────────────────────────┘
┌───────────────────────────┐
│    UNGROUPED_AGGREGATE    │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│        count_star()       │
└─────────────┬─────────────┘                             
┌─────────────┴─────────────┐
│         SEQ_SCAN          │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│             t             │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│   ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─   │
│           EC: 0           │
└───────────────────────────┘                             
D select count(*) from t;
┌──────────────┐
│ count_star() │
│    int64     │
├──────────────┤
│            0 │
└──────────────┘

@github-actions github-actions bot removed sql SQL Planner logical-expr Logical plan and expressions core Core DataFusion crate labels Jul 15, 2024
@tshauck
Copy link
Contributor Author

tshauck commented Jul 15, 2024

@jayzhan211 @alamb I've reverted this PR its original form. It adds a logical rule and updates the COUNT() function as a basis for discussion.

I also explored a little bit the idea of having a specific count_star function. The implementation is pretty simple and it's clear what's happening, though the biggest issue is it requires updating the Accumulator trait so that it can handle 0 param udafs. Or at least I don't think that's currently possible.

@jayzhan211
Copy link
Contributor

@tshauck Is it better to merge the logic of count-wild and count empty into single rule, given the similarity of them?

@alamb
Copy link
Contributor

alamb commented Jul 16, 2024

I will try and find time to review this PR tomorrow

@tshauck
Copy link
Contributor Author

tshauck commented Jul 17, 2024 via email

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @tshauck -- I think this looks good to me. I would prefer that we consolidate the optimizer passes, but we could also do it as part of a follow on PR (or never)

datafusion/functions-aggregate/src/count.rs Show resolved Hide resolved

/// Rewrite `Count()` to `Count(Expr:Literal(1))`.
#[derive(Default)]
pub struct CountEmptyRule {}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As each optimizer pass does have non trivial overhead (like it walks the plan trees) I think it would be better if this was combined into the existing CountWildcardRule if possible

@@ -103,10 +103,6 @@ SELECT power(1, 2, 3);
# Wrong window/aggregate function signature
#

# AggregateFunction with wrong number of arguments
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried to come up with some other query that has an invalid arguments for count(*) but I could not

@jayzhan211
Copy link
Contributor

As each optimizer pass does have non trivial overhead (like it walks the plan trees) I think it would be better if this was combined into the existing CountWildcardRule if possible

+1 on this, they are possible to be rewritten in one rule, it is ok to do it in another PR, but I hope we have only on rule at the end.

@tshauck
Copy link
Contributor Author

tshauck commented Jul 17, 2024

Thanks for the feedback @alamb @jayzhan211. I was unaware there was tangible overhead for each rule, so certainly makes sense to me to combine then. Still waiting on windows tests, but otherwise I've made the requested updates.

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Love it -- thank you @tshauck

@alamb alamb merged commit 0021356 into apache:main Jul 17, 2024
23 checks passed
@alamb
Copy link
Contributor

alamb commented Jul 17, 2024

Thanks for your guidance and encouragement @jayzhan211

Lordworms pushed a commit to Lordworms/arrow-datafusion that referenced this pull request Jul 23, 2024
* feat: add count empty rewrite

* feat: make count support zero args

* docs: add apache license

* tests: make count() valid

* tests: more tests

* refactor: sketch `AggregateFunctionPlanner`

* refactor: cleanup `AggregateFunctionPlanner`

* feat: add back rule

* Revert "feat: add back rule"

This reverts commit 2c4fc0a.

* Revert "refactor: cleanup `AggregateFunctionPlanner`"

This reverts commit 4550dbd.

* Revert "refactor: sketch `AggregateFunctionPlanner`"

This reverts commit 658671e.

* Apply suggestions from code review

Co-authored-by: Andrew Lamb <[email protected]>

* refactor: PR feedback

* style: fix indent

---------

Co-authored-by: Andrew Lamb <[email protected]>
wiedld pushed a commit to influxdata/arrow-datafusion that referenced this pull request Jul 31, 2024
* feat: add count empty rewrite

* feat: make count support zero args

* docs: add apache license

* tests: make count() valid

* tests: more tests

* refactor: sketch `AggregateFunctionPlanner`

* refactor: cleanup `AggregateFunctionPlanner`

* feat: add back rule

* Revert "feat: add back rule"

This reverts commit 2c4fc0a.

* Revert "refactor: cleanup `AggregateFunctionPlanner`"

This reverts commit 4550dbd.

* Revert "refactor: sketch `AggregateFunctionPlanner`"

This reverts commit 658671e.

* Apply suggestions from code review

Co-authored-by: Andrew Lamb <[email protected]>

* refactor: PR feedback

* style: fix indent

---------

Co-authored-by: Andrew Lamb <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Support COUNT() in addition to COUNT(*)
3 participants